-
Notifications
You must be signed in to change notification settings - Fork 1.1k
/
AbstractKafkaAvroDeserializer.java
605 lines (549 loc) · 23.1 KB
/
AbstractKafkaAvroDeserializer.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
/*
* Copyright 2018 Confluent Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.confluent.kafka.serializers;
import static io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils.getReflectData;
import static io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils.getReflectDataAllowNull;
import com.fasterxml.jackson.databind.JsonNode;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.CacheLoader;
import com.google.common.cache.LoadingCache;
import io.confluent.kafka.schemaregistry.client.rest.entities.RuleMode;
import java.io.InterruptedIOException;
import java.util.Collections;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.ExecutionException;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Type;
import org.apache.avro.generic.GenericContainer;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.specific.SpecificData;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificRecord;
import org.apache.kafka.common.errors.InvalidConfigurationException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.avro.reflect.ReflectData;
import org.apache.avro.reflect.ReflectDatumReader;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import io.confluent.kafka.schemaregistry.avro.AvroSchema;
import io.confluent.kafka.schemaregistry.avro.AvroSchemaProvider;
import io.confluent.kafka.schemaregistry.avro.AvroSchemaUtils;
import io.confluent.kafka.schemaregistry.client.rest.exceptions.RestClientException;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.header.Headers;
public abstract class AbstractKafkaAvroDeserializer extends AbstractKafkaSchemaSerDe {
private final DecoderFactory decoderFactory = DecoderFactory.get();
protected boolean useSpecificAvroReader = false;
protected Schema specificAvroReaderSchema = null;
protected boolean avroReflectionAllowNull = false;
protected boolean avroUseLogicalTypeConverters = false;
private final Map<String, Schema> readerSchemaCache = new ConcurrentHashMap<>();
private final LoadingCache<IdentityPair<Schema, Schema>, DatumReader<?>> datumReaderCache;
public AbstractKafkaAvroDeserializer() {
CacheLoader<IdentityPair<Schema, Schema>, DatumReader<?>> cacheLoader =
new CacheLoader<IdentityPair<Schema, Schema>, DatumReader<?>>() {
@Override
public DatumReader<?> load(IdentityPair<Schema, Schema> key) {
Schema writerSchema = key.getKey();
Schema readerSchema = key.getValue();
Schema finalReaderSchema = getReaderSchema(writerSchema, readerSchema);
boolean writerSchemaIsPrimitive =
AvroSchemaUtils.getPrimitiveSchemas().containsValue(writerSchema);
if (writerSchemaIsPrimitive) {
return new GenericDatumReader<>(writerSchema, finalReaderSchema,
avroUseLogicalTypeConverters
? AvroSchemaUtils.getGenericData()
: GenericData.get());
} else if (useSchemaReflection) {
return new ReflectDatumReader<>(writerSchema, finalReaderSchema,
avroUseLogicalTypeConverters
? getReflectData()
: ReflectData.get());
} else if (useSpecificAvroReader) {
return new SpecificDatumReader<>(writerSchema, finalReaderSchema);
} else {
return new GenericDatumReader<>(writerSchema, finalReaderSchema,
avroUseLogicalTypeConverters
? AvroSchemaUtils.getGenericData()
: GenericData.get());
}
}
};
datumReaderCache = CacheBuilder.newBuilder()
.maximumSize(DEFAULT_CACHE_CAPACITY)
.build(cacheLoader);
}
protected void configure(KafkaAvroDeserializerConfig config) {
configure(config, null);
}
/**
* Sets properties for this deserializer without overriding the schema registry client itself.
* Useful for testing, where a mock client is injected.
*/
protected void configure(KafkaAvroDeserializerConfig config, Class<?> type) {
configureClientProperties(config, new AvroSchemaProvider());
useSpecificAvroReader = config
.getBoolean(KafkaAvroDeserializerConfig.SPECIFIC_AVRO_READER_CONFIG);
if (useSpecificAvroReader && type != null) {
try {
specificAvroReaderSchema = ((SpecificRecord)type.getDeclaredConstructor().newInstance())
.getSchema();
} catch (Exception e) {
throw new ConfigException(
String.format(
"Error getting specificAvroReaderSchema from '%s'",
type.getName()
),
e
);
}
}
avroReflectionAllowNull = config
.getBoolean(KafkaAvroDeserializerConfig.AVRO_REFLECTION_ALLOW_NULL_CONFIG);
avroUseLogicalTypeConverters = config
.getBoolean(KafkaAvroSerializerConfig.AVRO_USE_LOGICAL_TYPE_CONVERTERS_CONFIG);
}
protected KafkaAvroDeserializerConfig deserializerConfig(Map<String, ?> props) {
return new KafkaAvroDeserializerConfig(props);
}
protected KafkaAvroDeserializerConfig deserializerConfig(Properties props) {
return new KafkaAvroDeserializerConfig(props);
}
/**
* Deserializes the payload without including schema information for primitive types, maps, and
* arrays. Just the resulting deserialized object is returned.
*
* <p>This behavior is the norm for Decoders/Deserializers.
*
* @param payload serialized data
* @return the deserialized object
*/
protected Object deserialize(byte[] payload) throws SerializationException {
return deserialize(null, isKey, payload, specificAvroReaderSchema);
}
/**
* Just like single-parameter version but accepts an Avro schema to use for reading
*
* @param payload serialized data
* @param readerSchema schema to use for Avro read (optional, enables Avro projection)
* @return the deserialized object
*/
protected Object deserialize(byte[] payload, Schema readerSchema) throws SerializationException {
return deserialize(null, isKey, payload, readerSchema);
}
protected Object deserialize(String topic, Boolean isKey, byte[] payload, Schema readerSchema)
throws SerializationException {
return deserialize(topic, isKey, null, payload, readerSchema);
}
protected Object deserialize(String topic, Boolean isKey, Headers headers,
byte[] payload, Schema readerSchema) throws SerializationException {
if (schemaRegistry == null) {
throw new InvalidConfigurationException(
"SchemaRegistryClient not found. You need to configure the deserializer "
+ "or use deserializer constructor with SchemaRegistryClient.");
}
if (payload == null) {
return null;
}
DeserializationContext context = new DeserializationContext(topic, isKey, headers, payload);
return context.read(context.schemaFromRegistry(),
readerSchema != null ? new AvroSchema(readerSchema) : null);
}
private Integer schemaVersion(String topic,
boolean isKey,
int id,
String subject,
AvroSchema schema,
Object result) {
try {
Integer version;
if (isDeprecatedSubjectNameStrategy(isKey)) {
subject = getSubjectName(topic, isKey, result, schema);
}
AvroSchema subjectSchema = (AvroSchema) schemaRegistry.getSchemaBySubjectAndId(subject, id);
version = schemaRegistry.getVersion(subject, subjectSchema);
return version;
} catch (InterruptedIOException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema version for id "
+ id;
throw new TimeoutException(errorMessage, e);
} catch (IOException e) {
throw new SerializationException("Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema version for id "
+ id, e);
} catch (RestClientException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema version for id "
+ id;
throw toKafkaException(e, errorMessage);
}
}
private String subjectName(String topic, boolean isKey, AvroSchema schemaFromRegistry) {
return isDeprecatedSubjectNameStrategy(isKey)
? null
: getSubjectName(topic, isKey, null, schemaFromRegistry);
}
/**
* Deserializes the payload and includes schema information, with version information from the
* schema registry embedded in the schema.
*
* @param payload the serialized data
* @return a GenericContainer with the schema and data, either as a {@link NonRecordContainer},
* {@link org.apache.avro.generic.GenericRecord}, or {@link SpecificRecord}
*/
protected GenericContainerWithVersion deserializeWithSchemaAndVersion(
String topic, boolean isKey, byte[] payload)
throws SerializationException, InvalidConfigurationException {
return deserializeWithSchemaAndVersion(topic, isKey, null, payload);
}
protected GenericContainerWithVersion deserializeWithSchemaAndVersion(
String topic, boolean isKey, Headers headers, byte[] payload)
throws SerializationException, InvalidConfigurationException {
// Even if the caller requests schema & version, if the payload is null we cannot include it.
// The caller must handle this case.
if (payload == null) {
return null;
}
// Annotate the schema with the version. Note that we only do this if the schema +
// version are requested, i.e. in Kafka Connect converters. This is critical because that
// code *will not* rely on exact schema equality. Regular deserializers *must not* include
// this information because it would return schemas which are not equivalent.
//
// Note, however, that we also do not fill in the connect.version field. This allows the
// Converter to let a version provided by a Kafka Connect source take priority over the
// schema registry's ordering (which is implicit by auto-registration time rather than
// explicit from the Connector).
DeserializationContext context = new DeserializationContext(topic, isKey, headers, payload);
AvroSchema schema = context.schemaForDeserialize();
Object result = context.read(schema,
specificAvroReaderSchema != null ? new AvroSchema(specificAvroReaderSchema) : null);
Integer version = schemaVersion(topic, isKey, context.getSchemaId(),
context.getSubject(), schema, result);
if (schema.rawSchema().getType().equals(Schema.Type.RECORD)) {
return new GenericContainerWithVersion((GenericContainer) result, version);
} else {
return new GenericContainerWithVersion(new NonRecordContainer(schema.rawSchema(), result),
version);
}
}
protected DatumReader<?> getDatumReader(Schema writerSchema, Schema readerSchema)
throws ExecutionException {
return datumReaderCache.get(new IdentityPair<>(writerSchema, readerSchema));
}
/**
* Normalizes the reader schema, puts the resolved schema into the cache.
* <li>
* <ul>if the reader schema is provided, use the provided one</ul>
* <ul>if the reader schema is cached for the writer schema full name, use the cached value</ul>
* <ul>if the writer schema is primitive, use the writer one</ul>
* <ul>if schema reflection is used, generate one from the class referred by writer schema</ul>
* <ul>if generated classes are used, query the class referred by writer schema</ul>
* <ul>otherwise use the writer schema</ul>
* </li>
*/
private Schema getReaderSchema(Schema writerSchema, Schema readerSchema) {
if (readerSchema != null) {
return readerSchema;
}
final boolean shouldSkipReaderSchemaCacheUsage = shouldSkipReaderSchemaCacheUsage(writerSchema);
if (!shouldSkipReaderSchemaCacheUsage) {
readerSchema = readerSchemaCache.get(writerSchema.getFullName());
}
if (readerSchema != null) {
return readerSchema;
}
boolean writerSchemaIsPrimitive =
AvroSchemaUtils.getPrimitiveSchemas().containsValue(writerSchema);
if (writerSchemaIsPrimitive) {
readerSchema = writerSchema;
} else if (useSchemaReflection) {
readerSchema = getReflectionReaderSchema(writerSchema);
readerSchemaCache.put(writerSchema.getFullName(), readerSchema);
} else if (useSpecificAvroReader) {
readerSchema = getSpecificReaderSchema(writerSchema);
if (!shouldSkipReaderSchemaCacheUsage) {
readerSchemaCache.put(writerSchema.getFullName(), readerSchema);
}
} else {
readerSchema = writerSchema;
}
return readerSchema;
}
private boolean shouldSkipReaderSchemaCacheUsage(Schema schema) {
return useSpecificAvroReader
&& (
schema.getType() == Type.ARRAY
|| schema.getType() == Type.MAP
|| schema.getType() == Type.UNION
);
}
@SuppressWarnings("unchecked")
private Schema getSpecificReaderSchema(Schema writerSchema) {
if (writerSchema.getType() == Type.ARRAY
|| writerSchema.getType() == Type.MAP
|| writerSchema.getType() == Type.UNION) {
return writerSchema;
}
Class<SpecificRecord> readerClass = SpecificData.get().getClass(writerSchema);
if (readerClass == null) {
throw new SerializationException("Could not find class "
+ writerSchema.getFullName()
+ " specified in writer's schema whilst finding reader's "
+ "schema for a SpecificRecord.");
}
try {
return readerClass.newInstance().getSchema();
} catch (InstantiationException e) {
throw new SerializationException(writerSchema.getFullName()
+ " specified by the "
+ "writers schema could not be instantiated to "
+ "find the readers schema.");
} catch (IllegalAccessException e) {
throw new SerializationException(writerSchema.getFullName()
+ " specified by the "
+ "writers schema is not allowed to be instantiated "
+ "to find the readers schema.");
}
}
private Schema getReflectionReaderSchema(Schema writerSchema) {
ReflectData reflectData = avroReflectionAllowNull
? (avroUseLogicalTypeConverters ? getReflectDataAllowNull() : ReflectData.AllowNull.get())
: (avroUseLogicalTypeConverters ? getReflectData() : ReflectData.get());
Class<?> readerClass = reflectData.getClass(writerSchema);
if (readerClass == null) {
throw new SerializationException("Could not find class "
+ writerSchema.getFullName()
+ " specified in writer's schema whilst finding reader's "
+ "schema for a reflected class.");
}
return reflectData.getSchema(readerClass);
}
class DeserializationContext {
private final String topic;
private final Boolean isKey;
private final Headers headers;
private final byte[] payload;
private final ByteBuffer buffer;
private final int schemaId;
DeserializationContext(
final String topic, final Boolean isKey, Headers headers, final byte[] payload) {
this.topic = topic;
this.isKey = isKey;
this.headers = headers;
this.payload = payload;
this.buffer = getByteBuffer(payload);
this.schemaId = buffer.getInt();
}
AvroSchema schemaFromRegistry() {
try {
String subjectName = isKey == null || strategyUsesSchema(isKey)
? getContext() : getSubject();
return (AvroSchema) schemaRegistry.getSchemaBySubjectAndId(subjectName, schemaId);
} catch (InterruptedIOException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId;
throw new TimeoutException(errorMessage, e);
} catch (IOException e) {
throw new SerializationException("Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId, e);
} catch (RestClientException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId;
throw toKafkaException(e, errorMessage);
}
}
AvroSchema schemaForDeserialize() {
try {
return isDeprecatedSubjectNameStrategy(isKey)
? AvroSchemaUtils.copyOf(schemaFromRegistry())
: (AvroSchema) schemaRegistry.getSchemaBySubjectAndId(getSubject(), schemaId);
} catch (InterruptedIOException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId;
throw new TimeoutException(errorMessage, e);
} catch (IOException e) {
throw new SerializationException("Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId, e);
} catch (RestClientException e) {
String errorMessage = "Error retrieving Avro "
+ getSchemaType(isKey)
+ " schema for id "
+ schemaId;
throw toKafkaException(e, errorMessage);
}
}
String getSubject() {
boolean usesSchema = strategyUsesSchema(isKey);
return subjectName(topic, isKey, usesSchema ? schemaFromRegistry() : null);
}
String getContext() {
return getContextName(topic);
}
String getTopic() {
return topic;
}
boolean isKey() {
return isKey;
}
int getSchemaId() {
return schemaId;
}
Object read(AvroSchema writerAvroSchema) {
return read(writerAvroSchema,
specificAvroReaderSchema != null ? new AvroSchema(specificAvroReaderSchema) : null);
}
Object read(AvroSchema writerAvroSchema, AvroSchema readerAvroSchema) {
try {
List<Migration> migrations = Collections.emptyList();
if (readerAvroSchema == null) {
if (metadata != null) {
readerAvroSchema = (AvroSchema) getLatestWithMetadata(getSubject());
} else if (useLatestVersion) {
readerAvroSchema =
(AvroSchema) lookupLatestVersion(getSubject(), writerAvroSchema, false);
}
if (readerAvroSchema != null) {
// set version on the writer schema
writerAvroSchema = schemaForDeserialize();
Integer version = schemaVersion(
topic, isKey, schemaId, getSubject(), writerAvroSchema, null);
writerAvroSchema = writerAvroSchema.copy(version);
migrations = getMigrations(getSubject(), writerAvroSchema, readerAvroSchema);
}
}
Schema writerSchema = writerAvroSchema.rawSchema();
Schema readerSchema = readerAvroSchema != null ? readerAvroSchema.rawSchema() : null;
DatumReader<?> reader;
if (!migrations.isEmpty()) {
// if migration is required, then initially use GenericDatumReader
reader = new GenericDatumReader<>(writerSchema, writerSchema,
avroUseLogicalTypeConverters
? AvroSchemaUtils.getGenericData()
: GenericData.get());
} else {
reader = getDatumReader(writerSchema, readerSchema);
}
int length = buffer.limit() - 1 - idSize;
Object result;
if (writerSchema.getType().equals(Schema.Type.BYTES)) {
byte[] bytes = new byte[length];
buffer.get(bytes, 0, length);
result = bytes;
} else {
int start = buffer.position() + buffer.arrayOffset();
result = reader.read(null, decoderFactory.binaryDecoder(buffer.array(),
start, length, null));
if (writerSchema.getType().equals(Schema.Type.STRING)) {
result = result.toString();
}
}
// First apply migration rules
if (!migrations.isEmpty()) {
result = executeMigrations(migrations, getSubject(), topic, headers, result);
}
if (readerAvroSchema == null) {
readerAvroSchema = writerAvroSchema;
}
if (result instanceof JsonNode) {
reader = getDatumReader(readerAvroSchema.rawSchema(), readerAvroSchema.rawSchema());
result = AvroSchemaUtils.toObject(
(JsonNode) result, readerAvroSchema, (DatumReader<Object>) reader);
}
// Next apply domain rules
result = executeRules(
getSubject(), topic, headers, payload, RuleMode.READ, null, readerAvroSchema, result
);
return result;
} catch (ExecutionException ex) {
throw new SerializationException("Error deserializing Avro message for id "
+ schemaId, ex.getCause());
} catch (RestClientException | IOException | RuntimeException e) {
// avro deserialization may throw AvroRuntimeException, NullPointerException, etc
throw new SerializationException("Error deserializing Avro message for id "
+ schemaId, e);
} finally {
postOp(payload);
}
}
}
static class IdentityPair<K, V> {
private final K key;
private final V value;
public IdentityPair(K key, V value) {
this.key = key;
this.value = value;
}
public K getKey() {
return key;
}
public V getValue() {
return value;
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
IdentityPair<?, ?> pair = (IdentityPair<?, ?>) o;
// Only perform identity check
return key == pair.key && value == pair.value;
}
@Override
public int hashCode() {
return System.identityHashCode(key) + System.identityHashCode(value);
}
@Override
public String toString() {
return "IdentityPair{"
+ "key=" + key
+ ", value=" + value
+ '}';
}
}
private static String getSchemaType(Boolean isKey) {
if (isKey == null) {
return "unknown";
} else if (isKey) {
return "key";
} else {
return "value";
}
}
}